Skip to content

Conversation

@andrewor14
Copy link
Contributor

What changes were proposed in this pull request?

This patch implements the CREATE TABLE command using the SessionCatalog. Previously we handled only CTAS and CREATE TABLE ... USING. This requires us to refactor CatalogTable to accept various fields (e.g. bucket and skew columns) and pass them to Hive.

WIP: Note that I haven't verified whether this actually works yet! But I believe it does.

How was this patch tested?

Tests will come in a future commit.

Andrew Or added 4 commits April 8, 2016 14:41
We need to reconcile the differences between what's added here in
SparkSqlParser and HiveSqlParser. That will come in the next
commit.

This currently still fails tests, obviously because create table
is not implemented yet!
Before: CatalogTable has schema, partitionColumns and sortColumns.
There are no constraints between the 3. However, Hive will
complain if schema and partitionColumns overlap.

After: CatalogTable has schema, partitionColumnNames,
sortColumnNames, bucketColumnNames and skewColumnNames. All the
columns must be a subset of schema. This means splitting up
schema into (schema, partitionCols) before passing it to Hive.

This allows us to store the columns more uniformly. Otherwise
partition columns would be the odd one out. This commit also
fixes "alter table bucketing", which was incorrectly using
partition columns as bucket columns.
This involves reverting part of the changes in an earlier commit,
where we tried to implement the parsing logic in the general SQL
parser and introduced a bunch of case classes that we won't end
up using.

As of this commit the actual CREATE TABLE logic is not there yet.
It will come in a future commit.
* ALTER TABLE table1 RENAME TO table2;
* ALTER VIEW view1 RENAME TO view2;
* }}}
*/
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

moved to tables.scala. I just moved this one command for now to avoid inflating the diff too much.


fileFormat
: INPUTFORMAT inFmt=STRING OUTPUTFORMAT outFmt=STRING (SERDE serdeCls=STRING)?
(INPUTDRIVER inDriver=STRING OUTPUTDRIVER outDriver=STRING)? #tableFileFormat
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@hvanhovell I deleted the INPUTDRIVER and OUTPUTDRIVER here because Hive doesn't support it. Why was this added in the first place? Is there any supporting documentation for this somewhere?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@andrewor14 I wanted to make sure we supported the same grammar as Hive and I used their grammars as a basis. So this is defined in the following two locations:

The main idea was that I could throw better errors. But if it is not supported by Hive itself then please remove it!

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's remove it. I have never seen this before and it is not documented anywhere.

@SparkQA
Copy link

SparkQA commented Apr 9, 2016

Test build #55425 has finished for PR 12271 at commit 5e0fe03.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 9, 2016

Test build #55426 has finished for PR 12271 at commit f7501d9.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

@andrewor14 Just want to let you know, @xwu0226 is doing the command SHOW CREATE TABLE. He is writing many test cases in the PR: #12132. It might help you in this PR. Thanks!

* [COMMENT table_comment]
* [PARTITIONED BY (col3 data_type [COMMENT col_comment], ...)]
* [CLUSTERED BY (col1, ...) [SORTED BY (col1 [ASC|DESC], ...)] INTO num_buckets BUCKETS]
* [SKEWED BY (col1, col2, ...) ON ((col_value, col_value, ...), ...)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@andrewor14 I just did a quick check with our InsertIntoHiveTable command. Seems this command does not really understand how to handle a table having specifications on CLUSTERED BY, SORTED BY or SKEWED BY. How about we just throw exceptions when a define provide these specs?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

@andrewor14 andrewor14 changed the title [SPARK-14388][SQL][WIP] Implement CREATE TABLE [SPARK-14388][SQL] Implement CREATE TABLE Apr 11, 2016
@SparkQA
Copy link

SparkQA commented Apr 12, 2016

Test build #55552 has finished for PR 12271 at commit 2e95ecf.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@andrewor14
Copy link
Contributor Author

I'll have a look at the failing tests in a couple of hours.

* [COMMENT table_comment]
* [PARTITIONED BY (col3 data_type [COMMENT col_comment], ...)]
* [CLUSTERED BY (col1, ...) [SORTED BY (col1 [ASC|DESC], ...)] INTO num_buckets BUCKETS]
* [SKEWED BY (col1, col2, ...) ON ((col_value, col_value, ...), ...) [STORED AS DIRECTORIES]]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove this line while resolving the conflicts?

Andrew Or added 2 commits April 12, 2016 16:42
Previously we always converted the data type string to lower case.
However, for struct fields this also converts the struct field
names to lower case. This is not what tests (or perhaps user code)
expects.
@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55658 has finished for PR 12271 at commit 50a2054.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55666 has finished for PR 12271 at commit 8dc554a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@andrewor14
Copy link
Contributor Author

I've ignored some tests in HiveCompatibilitySuite for now. I'll have a look at them shortly. I just wanted to see if the rest of the tests will pass.

@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55674 has finished for PR 12271 at commit 8e273fd.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

"date_join1",
"date_serde",
"decimal_1",
//"decimal_1", // TODO: cannot parse column decimal(5)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea. We should support decimal(5). At here, the scale is 0.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

// just convert the whole type string to lower case, otherwise the struct field names
// will no longer be case sensitive. Instead, we rely on our parser to get the proper
// case before passing it to Hive.
HiveMetastoreTypes.toDataType(col.dataType.getText).simpleString,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we need to use parseDataType provided in AbstractSqlParser.

Andrew Or added 3 commits April 12, 2016 23:26
There were a few differences in DESCRIBE TABLE:
- output format should be HiveIgnoreKeyTextOutputFormat
- num buckets should be -1
- last access time should be -1
- EXTERNAL should not be set to false for managed table

After making these changes out result now matches Hive's.
CatalystSqlParser knows how to parse decimal(5)!
// just convert the whole type string to lower case, otherwise the struct field names
// will no longer be case sensitive. Instead, we rely on our parser to get the proper
// case before passing it to Hive.
CatalystSqlParser.parseDataType(col.dataType.getText).simpleString,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MINOR/NIT: The DataType parsing is done in the AstBuilder, so we really don't need to parse this string again. You could use (magical/evil) typedVisit[DataType](col.dataType) here.

@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55692 has finished for PR 12271 at commit 59edce3.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55693 has finished for PR 12271 at commit a60e66a.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55696 has finished for PR 12271 at commit 02738fe.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

table.storage.outputFormat.map(toOutputFormat).foreach(hiveTable.setOutputFormatClass)
table.storage.serde.foreach(hiveTable.setSerializationLib)
hiveTable.setSerializationLib(
table.storage.serde.getOrElse("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hmm... Without this PR, the storage related fields of the table defined in org.apache.spark.sql.hive.MetastoreDataSourcesSuite.persistent JSON table are:

[SerDe Library:         org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe    ]
[InputFormat:           org.apache.hadoop.mapred.SequenceFileInputFormat     ]
[OutputFormat:          org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat    ]

Looks like the change at here somehow breaks the test (without the change, we can somehow trigger a weird code path in Hive and get at least have one column called col).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For now, if the schema does not have any field, can we set it to the following one?

[# col_name             data_type               comment             ]
[        ]
[col                    array<string>           from deserializer   ]

So, we try to preserve the existing (and weird) behavior.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I submitted PR to try it out (#12363). You can find my change in the last commit (ab70cb7).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why are there all these undocumented implicit behaviors in Hive :(

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now, after a few bad experiences, I think we need to add all the test cases to ensure the Hive APIs work as what we expected. See another issue I hit in alter table ... drop partition: #12220 (diff)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I feel it is better to have more checks in SessionCatalog to make sure that a request is valid (it can help your case).

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, agree if you are talking about my case. : )

@andrewor14
Copy link
Contributor Author

OK, I cherry-picked your changes and updated the comment.

@SparkQA
Copy link

SparkQA commented Apr 13, 2016

Test build #55727 has finished for PR 12271 at commit 55957bd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yhuai
Copy link
Contributor

yhuai commented Apr 13, 2016

OK. Merging to master.

@asfgit asfgit closed this in 7d2ed8c Apr 13, 2016
@andrewor14 andrewor14 deleted the create-table-ddl branch June 22, 2016 17:48
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants